6c89c6
@@ -30,15 +30,6 @@
import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.lmax.disruptor.BlockingWaitStrategy;
-import com.lmax.disruptor.EventHandler;
-import com.lmax.disruptor.ExceptionHandler;
-import com.lmax.disruptor.LifecycleAware;
-import com.lmax.disruptor.TimeoutException;
-import com.lmax.disruptor.dsl.Disruptor;
-import com.lmax.disruptor.dsl.ProducerType;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -68,6 +59,15 @@
import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.ExceptionHandler;
+import com.lmax.disruptor.LifecycleAware;
+import com.lmax.disruptor.TimeoutException;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
 /**
  * The default implementation of FSWAL.
  */
@@ -499,6 +499,7 @@
public class FSHLog extends AbstractFSWAL<Writer> {
     private volatile long sequence;
     // Keep around last exception thrown. Clear on successful sync.
     private final BlockingQueue<SyncFuture> syncFutures;
+    private volatile SyncFuture takeSyncFuture = null;
 
     /**
      * UPDATE!
@@ -546,6 +547,7 @@
public class FSHLog extends AbstractFSWAL<Writer> {
       if (!syncFuture.done(currentSequence, t)) {
         throw new IllegalStateException();
       }
+
       // This function releases one sync future only.
       return 1;
     }
@@ -589,13 +591,21 @@
public class FSHLog extends AbstractFSWAL<Writer> {
       return sequence;
     }
 
+    boolean areSyncFuturesReleased() {
+      // check whether there is no sync futures offered, and no in-flight sync futures that is being
+      // processed.
+      return syncFutures.size() <= 0
+          && takeSyncFuture == null;
+    }
+
     public void run() {
       long currentSequence;
       while (!isInterrupted()) {
         int syncCount = 0;
-        SyncFuture takeSyncFuture;
+
         try {
           while (true) {
+            takeSyncFuture = null;
             // We have to process what we 'take' from the queue
             takeSyncFuture = this.syncFutures.take();
             currentSequence = this.sequence;
@@ -975,11 +985,23 @@
public class FSHLog extends AbstractFSWAL<Writer> {
      * @return True if outstanding sync futures still
      */
     private boolean isOutstandingSyncs() {
+      // Look at SyncFutures in the EventHandler
       for (int i = 0; i < this.syncFuturesCount; i++) {
         if (!this.syncFutures[i].isDone()) {
           return true;
         }
       }
+
+      return false;
+    }
+
+    private boolean isOutstandingSyncsFromRunners() {
+      // Look at SyncFutures in the SyncRunners
+      for (SyncRunner syncRunner: syncRunners) {
+        if(syncRunner.isAlive() && !syncRunner.areSyncFuturesReleased()) {
+          return true;
+        }
+      }
       return false;
     }
 
@@ -1095,11 +1117,13 @@
public class FSHLog extends AbstractFSWAL<Writer> {
         // Wait on outstanding syncers; wait for them to finish syncing (unless we've been
         // shutdown or unless our latch has been thrown because we have been aborted or unless
         // this WAL is broken and we can't get a sync/append to complete).
-        while (!this.shutdown && this.zigzagLatch.isCocked()
+        while ((!this.shutdown && this.zigzagLatch.isCocked()
             && highestSyncedTxid.get() < currentSequence &&
             // We could be in here and all syncs are failing or failed. Check for this. Otherwise
             // we'll just be stuck here for ever. In other words, ensure there syncs running.
-        isOutstandingSyncs()) {
+            isOutstandingSyncs())
+            // Wait for all SyncRunners to finish their work so that we can replace the writer
+            || isOutstandingSyncsFromRunners()) {
           synchronized (this.safePointWaiter) {
             this.safePointWaiter.wait(0, 1);
           }
